[fix][broker] Fix compacted read could be stuck forever or message loss due to cursor mark delete#25998
Merged
BewareMyPower merged 6 commits intoJun 12, 2026
Conversation
…ss due to delayed acknowledgment
poorbarcode
approved these changes
Jun 11, 2026
Contributor
Author
|
|
poorbarcode
reviewed
Jun 11, 2026
Contributor
Author
|
After removing |
Contributor
Author
|
Oh I've figured out the reason, it's because both system topic reader and the normal reader have |
codelipenghui
approved these changes
Jun 11, 2026
dao-jun
approved these changes
Jun 11, 2026
void-ptr974
reviewed
Jun 12, 2026
void-ptr974
approved these changes
Jun 12, 2026
void-ptr974
left a comment
Contributor
There was a problem hiding this comment.
Thanks for the fix! The updated change looks good to me.
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Motivation
We observed the system topic reader encountered a typical bug case:
hasMessageAvailable()returns true butreadNext()is not able to read any message. It causes all topics in the same namespace fail to load.After digging into details, I've figured out the cause for this incident. During the incident, there are two unload operations on bundles of the same namespace in short time:
__change_eventstopicbroker-0, the message id isA:2__change_eventstopic frombroker-0tobroker-1broker-1, withA:2as the start message id.Then the reader was stuck. From the heap dump, I found the following important info from the dispatcher:
As you can see, all entries are compacted (the original ledger id is
A), the managed ledger was just empty with an empty ledger whose id isB.From the logs, we can see the cursor's initial mark delete position and read position is different:
But the
ackPosisA:2in the heap dump, andreadPosisB:0. I also checked the topic compaction context's cache and dumped the compacted ledger to verify there is even nofindStartPointcall on the compacted ledger.The root cause is:
broker-0, it stored the message IDA:2toPersistentAcknowledgmentsGroupingTracker#lastCumulativeAckviaacknowledgeCumulativeAsync.acknowledgmentGroupTime), the reader had already switched the connection tobroker-1, then it flushed the ACK request tobroker-1broker-1received the ACK request and simply calledcursor.asyncMarkDelete(A:2).pulsar/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java
Line 474 in a045e4c
The mark delete position was simply set to
A:2:pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java
Line 2216 in a045e4c
However, the read position was advanced to the next valid position (
B:0) in the managed ledger:pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java
Line 2225 in a045e4c
while the compaction horizon is just
B:-1, which is behind the new read position:pulsar/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java
Lines 55 to 56 in a045e4c
and the read operation went through the managed cursor directly and would be stuck forever.
Modifications
When a subscription has a compacted consumer, if the acknowledged position's ledger id does not exist in the managed ledger, just do nothing. Otherwise, even if the next valid ledger has messages to read, the compacted entries could be skipped, which leads to message loss.
Verifying this change
testReceiveAckAfterReconnectionOnEmptyLedgercovers the case exactly.Others
Consumer does not have this issue, even if it's non-durable, because consumer's
startMessageIdisnull, so inasyncReadCompactedEntries, in the first read,readFromEarliestis always true.